155a64
@@ -2421,25 +2421,22 @@
public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
   /**
    * Perform a batch of mutations.
    * It supports only Put and Delete mutations and will ignore other types passed.
-   * @param mutations the list of mutations
+   * @param batchOp contains the list of mutations
    * @return an array of OperationStatus which internally contains the
    *         OperationStatusCode and the exceptionMessage if any.
    * @throws IOException
    */
   OperationStatus[] batchMutate(BatchOperationInProgress<?> batchOp) throws IOException {
     boolean initialized = false;
-    while (!batchOp.isDone()) {
-      if (!batchOp.isInReplay()) {
-        checkReadOnly();
-      }
-      checkResources();
-
-      long newSize;
-      Operation op = Operation.BATCH_MUTATE;
-      if (batchOp.isInReplay()) op = Operation.REPLAY_BATCH_MUTATE;
-      startRegionOperation(op);
+    Operation op = batchOp.isInReplay() ? Operation.REPLAY_BATCH_MUTATE : Operation.BATCH_MUTATE;
+    startRegionOperation(op);
+    try {
+      while (!batchOp.isDone()) {
+        if (!batchOp.isInReplay()) {
+          checkReadOnly();
+        }
+        checkResources();
 
-      try {
         if (!initialized) {
           this.writeRequestsCount.add(batchOp.operations.length);
           if (!batchOp.isInReplay()) {
@@ -2448,13 +2445,13 @@
public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
           initialized = true;
         }
         long addedSize = doMiniBatchMutation(batchOp);
-        newSize = this.addAndGetGlobalMemstoreSize(addedSize);
-      } finally {
-        closeRegionOperation(op);
-      }
-      if (isFlushSize(newSize)) {
-        requestFlush();
+        long newSize = this.addAndGetGlobalMemstoreSize(addedSize);
+        if (isFlushSize(newSize)) {
+          requestFlush();
+        }
       }
+    } finally {
+      closeRegionOperation(op);
     }
     return batchOp.retCodeDetails;
   }
@@ -2583,7 +2580,7 @@
public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
         boolean shouldBlock = numReadyToWrite == 0;
         RowLock rowLock = null;
         try {
-          rowLock = getRowLock(mutation.getRow(), shouldBlock);
+          rowLock = getRowLockInternal(mutation.getRow(), shouldBlock);
         } catch (IOException ioe) {
           LOG.warn("Failed getting lock in batch put, row="
             + Bytes.toStringBinary(mutation.getRow()), ioe);
@@ -3767,45 +3764,53 @@
public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
    * @throws IOException if waitForLock was true and the lock could not be acquired after waiting
    */
   public RowLock getRowLock(byte[] row, boolean waitForLock) throws IOException {
-    checkRow(row, "row lock");
     startRegionOperation();
     try {
-      HashedBytes rowKey = new HashedBytes(row);
-      RowLockContext rowLockContext = new RowLockContext(rowKey);
+      return getRowLockInternal(row, waitForLock);
+    } finally {
+      closeRegionOperation();
+    }
+  }
 
-      // loop until we acquire the row lock (unless !waitForLock)
-      while (true) {
-        RowLockContext existingContext = lockedRows.putIfAbsent(rowKey, rowLockContext);
-        if (existingContext == null) {
-          // Row is not already locked by any thread, use newly created context.
-          break;
-        } else if (existingContext.ownedByCurrentThread()) {
-          // Row is already locked by current thread, reuse existing context instead.
-          rowLockContext = existingContext;
-          break;
-        } else {
-          if (!waitForLock) {
-            return null;
-          }
-          try {
-            // Row is already locked by some other thread, give up or wait for it
-            if (!existingContext.latch.await(this.rowLockWaitDuration, TimeUnit.MILLISECONDS)) {
-              throw new IOException("Timed out waiting for lock for row: " + rowKey);
-            }
-          } catch (InterruptedException ie) {
-            LOG.warn("Thread interrupted waiting for lock on row: " + rowKey);
-            InterruptedIOException iie = new InterruptedIOException();
-            iie.initCause(ie);
-            throw iie;
+  /**
+   * A version of getRowLock(byte[], boolean) to use when a region operation has already been
+   * started (the calling thread has already acquired the region-close-guard lock).
+   */
+  protected RowLock getRowLockInternal(byte[] row, boolean waitForLock) throws IOException {
+    checkRow(row, "row lock");
+    HashedBytes rowKey = new HashedBytes(row);
+    RowLockContext rowLockContext = new RowLockContext(rowKey);
+
+    // loop until we acquire the row lock (unless !waitForLock)
+    while (true) {
+      RowLockContext existingContext = lockedRows.putIfAbsent(rowKey, rowLockContext);
+      if (existingContext == null) {
+        // Row is not already locked by any thread, use newly created context.
+        break;
+      } else if (existingContext.ownedByCurrentThread()) {
+        // Row is already locked by current thread, reuse existing context instead.
+        rowLockContext = existingContext;
+        break;
+      } else {
+        if (!waitForLock) {
+          return null;
+        }
+        try {
+          // Row is already locked by some other thread, give up or wait for it
+          if (!existingContext.latch.await(this.rowLockWaitDuration, TimeUnit.MILLISECONDS)) {
+            throw new IOException("Timed out waiting for lock for row: " + rowKey);
           }
+        } catch (InterruptedException ie) {
+          LOG.warn("Thread interrupted waiting for lock on row: " + rowKey);
+          InterruptedIOException iie = new InterruptedIOException();
+          iie.initCause(ie);
+          throw iie;
         }
       }
-
-      // allocate new lock for this thread
-      return rowLockContext.newLock();
-    } finally {
-      closeRegionOperation();
     }
+
+    // allocate new lock for this thread
+    return rowLockContext.newLock();
   }
 
   /**
